Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ZOOKEEPER-4541 Ephemeral znode owned by closed session visible in 1 of 3 servers #1925

Closed
wants to merge 16 commits into from

Conversation

jonmv
Copy link
Contributor

@jonmv jonmv commented Sep 23, 2022

This fixes two bugs in shutdown logic, in the zookeeper server.

  1. The SendAckRequestProcessor may die when attempting to close its Learner owner's socket, to signal that something went wrong, if the learner already closed the socket because something (the same thing) went wrong (namely, the leader disconnecting). This is fixed by simply checking for nullity.
  2. ZooKeeperServer.shutdown(boolean) is not present in child classes, so many uses here fail to properly shut down child resources, such as the SyncRequestProcessor. This is fixed by refactoring shutdown for the child classes.

A unit test is also added, that fails when either of the two fixes are not present.
To be precise, it fails only because the SyncRequestProcessor is never shut down (thread leak), once the first fix is applied; I didn't spend more time looking for other weird failures that may arise from what is obviously a bug anyway.

See ZOOKEEPER-4541 for full details.

@eolivelli eolivelli changed the title Jonmv/zookeeper 4541 ZOOKEEPER-4541 Ephemeral znode owned by closed session visible in 1 of 3 servers Sep 23, 2022
@jonmv
Copy link
Contributor Author

jonmv commented Sep 23, 2022

@hanm I believe you review a related PR earlier, so perhaps you're the right reviewer here as well?

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

Great work.
Did you experiment this fix in some test environment ?

@eolivelli
Copy link
Contributor

I don't know why I cannot add @hanm as "reviewer", btw I hope he will receive the notification

@@ -64,7 +66,8 @@ public void flush() throws IOException {
} catch (IOException e) {
LOG.warn("Closing connection to leader, exception during packet send", e);
try {
if (!learner.sock.isClosed()) {
Socket socket = learner.sock;
if ( socket != null && ! learner.sock.isClosed()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should probably use socket in the second condition too? In case it changes after the first check.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, that was of course the intention :) Fixed!

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hi jonmv, i have read the jira-4541. I am confusing. ZK1 does not send ack to leader , ZK1 recieves commit from leader. It seems not conform ZAB protocol. Please help me figure out, thanks

@jonmv
Copy link
Contributor Author

jonmv commented Sep 23, 2022

LGTM

Great work. Did you experiment this fix in some test environment ?

Not yet, but we may do that before we merge, if you wish. We will find a way to run a patched 3.8.0, probably early next week.

@jonmv
Copy link
Contributor Author

jonmv commented Sep 23, 2022

Thanks for the quick reply!

@jonmv
Copy link
Contributor Author

jonmv commented Sep 30, 2022

We've had this running for almost a week now, without any issues, and the data inconsistencies have not been observed. The sample size isn't large enough to conclude, though :)

Anyway, we saw some other digest mismatches, and I started digging around for their cause. I found one problem introduced with this commit, fixed in 8121711.
The problem was a COMMIT between NEWLEADER (which flushes the packetsNotCommited) and UPTODATE would crash the learner, which would peek at this queue and expect entries in it. This is fixed by not clearing the packetsNotCommited on NEWLEADER; instead, the already written entries are simply skipped when updating the log after UPTODATE.

Working on the above, I also found the fix for reconfig between NEWLEADER and UPTODATE, in this commit, to be incomplete: since the packetsNotCommited is no longer emptied after NEWLEADER, the head doesn't change, and if there are other PROPOSALs between the NEWLEADER and the PROPOSAL that the COMMITANDACTIVATE is meant for, then reconfig still doesn't happen. The unit test added back then was insufficient to test this. My fix to this is in f24eb51, and is to simply traverse the packetsNotCommitted, looking for a matching Zxid. I can't imaging this being a performance issue.

@sonatype-lift
Copy link

sonatype-lift bot commented Sep 30, 2022

⚠️ 52 God Classes were detected by Lift in this project. Visit the Lift web console for more details.

@jonmv
Copy link
Contributor Author

jonmv commented Sep 30, 2022

The symptom of the bug fixed in the first of these two commits is that the learner crashes with an NPE during sync, and then, when it restarts, it typically writes a duplicated series of transactions to its transaction log, complains about that, and later may observe a digest mismatch when replaying that transaction log from file, during startup.

@jeffrey-xiao
Copy link

jeffrey-xiao commented Sep 30, 2022

Ah, I think we raced -- I just minted a PR to resolve ZOOKEEPER-4394 which is the problem you described. I'm happy with either solution, but perhaps you can also take a look at my PR?

@jonmv
Copy link
Contributor Author

jonmv commented Oct 1, 2022

We raced indeed :) Given that this PR addresses some additional concerns, I'd vote for it to be merged.

@jonmv
Copy link
Contributor Author

jonmv commented Oct 3, 2022

Actually, your version is better. I'm incorporating that here instead, if you don't mind? I still think it's a good idea to ensure the pending writes are actually flushed, before ack'ing the NEWLEADER, both because we should ensure they're on persistent storage before ack'ing, and also because that reduces the otherwise random order of ACKs the leader would observe, when SyncRequestProcessor.run races with Learner.syncWithLeader.

@jonmv jonmv force-pushed the jonmv/ZOOKEEPER-4541 branch 3 times, most recently from abaf8af to 4aab1fd Compare October 3, 2022 10:35
@jonmv
Copy link
Contributor Author

jonmv commented Oct 3, 2022

Hmm, no, this isn't quite right either, although none of the tests fail.
Not sure why the rendezvous-with-sync-thread didn't work, and that's probably the right way to do this. It could be just insufficient test-setup, of course.

@jonmv
Copy link
Contributor Author

jonmv commented Oct 3, 2022

Hmm, no, the code has a race, as it is now. The LeaderHandler expects the first ACK after starting a DIFF sync to be the NEWLEADER ack, but if there are lots of PROPOSALs in the diff, before the NEWLEADER, then this may also cause an ACK to be sent, which will crash the Leader.wantForNewLeaderAck.

Ensuring the transactions are indeed flushed (through the usual request processor pipeline) guarantees these ACKs, and thus always crashes the leader. Meh.

@jonmv
Copy link
Contributor Author

jonmv commented Oct 3, 2022

There ... Not pretty, but seems like the only way to make this right is by making it possible to delay the ACKs that would otherwise be sent once the SyncRequestProcessor flushes, so they arrive after the ACK og the NEWLEADER.
This is implemented by adding additional special requests that can be enqueued by the sync processor—one to rendezvous with it, from the syncing thread at startup, to ensure TXNs are actually flushed; and two to toggle whether to delay forwarding to the SendAckRequestProcessor, used by the FollowerZooKeeperServer.

@jeffrey-xiao
Copy link

FWIW, your recent commit looks good to me.

Another possible approach is to bypass the request processor pipeline entirely like what was done in #1848.

if (zk instanceof FollowerZooKeeperServer) {
FollowerZooKeeperServer fzk = (FollowerZooKeeperServer) zk;
fzk.syncProcessor.setDelayForwarding(false);
fzk.syncProcessor.syncFlush();

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the reason why we need the second syncFlush?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To ensure consistent ordering of the UPTODATE ACK, vs ACKs from PROPOSALs. The real leader doesn't care, but unit tests may.

@jonmv
Copy link
Contributor Author

jonmv commented Oct 19, 2022

So ... you may be right we don't need to keep all this auxiliary structure during sync, but I believe we need to if we want to precisely preserve today's behaviour (except what's needed to fixi those bugs, obviously).
What complicates the sync is that some transactions aren't logged and ack'ed; and possibly also that state is used while syncing. I can't say whether the first behaviour is needed, and whether the second is actually the case, without a much deeper dive into all of this, but I do see tests failing when I change this behaviour.
Perhaps it is a good idea to first fix these bugs, and then possibly look for ways to simplify?

@jonmv
Copy link
Contributor Author

jonmv commented Jan 3, 2023

Any further thoughts on this @breed, @eolivelli ?

@jonmv
Copy link
Contributor Author

jonmv commented Jan 23, 2023

I could add that after patching with these commits (not the last one, which should be purely refactoring), we've had zero issues with inconsistent ZK clusters. This is across several hundred thousand rolling cluster restarts. Previously, we typically had one or two broken clusters each week, and had to intervene manually in each case, i.e., a one-in-thousand chance of breaking across a restart.

@jeffrey-xiao
Copy link

As another data point, we're also running multiple clusters with weekly restarts and have not seen issues with inconsistent ZK clusters with this patch. I am very interested in getting this merged and was wondering what's left to push this PR through?

IMO, this is pretty high priority bug to fix because it requires manual intervention to recover from. Otherwise, the cluster is in a permanently inconsistent state.

Copy link

@jeffrey-xiao jeffrey-xiao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As a meta note, I wonder if it's worth splitting this PR into multiple PRs because it fixes distinct bugs (ZOOKEEPER-4409, ZOOKEEPER-4502, ZOOKEEPER-4394, ZOOKEEPER-4541). Likely not worth the effort, but perhaps it will get through review more easily that way ;)

@jonmv
Copy link
Contributor Author

jonmv commented Feb 3, 2023

As a meta note, I wonder if it's worth splitting this PR into multiple PRs because it fixes distinct bugs (ZOOKEEPER-4409, ZOOKEEPER-4502, ZOOKEEPER-4394, ZOOKEEPER-4541). Likely not worth the effort, but perhaps it will get through review more easily that way ;)

Oof, would probably have been a good idea, but one thing led to another, and now it'd be a lot of work to split this 😬 🙂

@jonmv
Copy link
Contributor Author

jonmv commented Mar 16, 2023

@eolivelli any thoughts on what to do next here? I think it would be good to conclude this work soon. It's getting a bit stale :)

Copy link
Contributor

@fanyang89 fanyang89 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi, jonmv. I'm very interested in your work and read it carefully.
Some questions are coming below, maybe we could discuss them.

@@ -106,6 +106,8 @@ protected void setupRequestProcessors() {
if (syncRequestProcessorEnabled) {
syncProcessor = new SyncRequestProcessor(this, null);
syncProcessor.start();
} else {
syncProcessor = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

syncProcessor as an ObserverZooKeeperServer field should have a default value of null.
Does setting null here makes a difference?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, I'm just used to always assigning (to final fields). This can be removed again.

@@ -174,6 +224,21 @@ public void run() {
break;
}

if (si == turnForwardingDelayOn) {
nextProcessor.close();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At the ctor of SyncRequestProcessor, nextProcessor may be null.
Can this be an NPE at ObserverZooKeeperServer(with syncRequestProcessorEnabled=true)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Only followers enqueue these special requests, so that can't happen. Observers don't ack txns, as far as I remember?

continue;
}
if (si == turnForwardingDelayOff) {
nextProcessor.open();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This naming here is confusing.
The intention here is: to get the turning delaying off request, open the gate, then flush all pending requests to the downstream processor.
nextProcessor.open() is to open the gate or turn the delay on?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

open() opens the gate. What about startDelaying() and flushAndStopDelaying()?

while (createZxid1 != follower.fzk.getLastProcessedZxid() && System.currentTimeMillis() < doom) {
Thread.sleep(1);
}
assertEquals(createZxid1, follower.fzk.getLastProcessedZxid());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Running the unit test shows that this assertion is not always true.
After txn(1, counter=3) is flushed, SyncRequestProcessor can take() txn(1, 4) and add to the toFlush queue without a flush(), then poll() returns a null, and the processor flushes. Txn(1,4) may or may not flush; it depends on the order.
A simple workaround is to enable flushDelay (via zookeeper.flushDelay) so that a flush for txn(1,4) is not called from an incoming null request. Add a barrier maybe?
It's likely to happen in an earlier version of the JDK(e.g., 1.8, 10, 11, etc., it has yet to occur at JDK18 in my test env, but why?)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, thanks for spotting this. The intention was to wait for the second create txn ID, not the first.

The read variable is volatile, so changes should be visible, given enough (sleep) time to let the other thread do its work.
The construct was copied from a different test in the same class (lines 1101–1107), so if this is still unstable, I'd expect that test to also need an update.

@kezhuw
Copy link
Member

kezhuw commented May 4, 2023

Hmm, that other PR doesn't look right to me. It fails to store the TXNs that aren't already committed before ACKing the NEWLEADER, which was what ZOOKEEPER-3911 was all about, in the first place. Agree?

// ZOOKEEPER-3911: make sure sync the uncommitted logs before commit them (ACK NEWLEADER).

Given that, I would say #1445 itself is misleading or we are misunderstanding the jira part of ZOOKEEPER-3911, or both. It should sync only txns in DIFF sync, all proposals in this phase are considered committed by new leader. I think ZOOKEEPER-4394 already/almost listed the point: NEWLEADER is not appended intermediately/atomically after proposals in DIFF sync. I guess we are not aware of this by the time of #1445.

These ongoing(committed or not) proposals are simply beyond of discussion, these belongs to broadcast phase(See Zab, Zab1.0). I considered them as gap between paper and implementation. Steps in paper are mostly like atomic, but implementations are not.

Back to this pr, the good part is that it is verified in production. The bad part is that it is giant and mixed several issues and areas.

Personally, I would suggest to fix above issues separately. There are other possible issues in synchronization phase.

  • ZOOKEEPER-4643: Committed txns may be improperly truncated if follower crashes right after updating currentEpoch but before persisting txns to disk
  • NEWLEADER is not sent intermediately after DIFF syncs. This diverges from the paper. ZOOKEEPER-4394 pointed out this.
  • It is not guaranteed/asserted in LearnerHandler that the first ACK is targeting NEWLEADER. ZOOKEEPER-4685 pointed out this.
  • Unmatched commits are causing log.warns in synchronization phase, while they cause system exit in FollowerZooKeeperServer.commit.

IMO, this is pretty high priority bug to fix because it requires manual intervention to recover from. Otherwise, the cluster is in a permanently inconsistent state.

I think it would be good to conclude this work soon. It's getting a bit stale :)

Hmm, some of my prs #1820(merged), #1859(approved) and #1847 are all almost one year from its creation. I also opened apache/bookkeeper#3041(data loss, 1 year), apache/pulsar#7490(data duplication, 3 years) and google/leveldb#375(data inconsistency, 3 years until google/leveldb#339 got merged). I guess we should believe in time 😮‍💨.

Oof, would probably have been a good idea, but one thing led to another, and now it'd be a lot of work to split this 😬 🙂

Maybe we can start from fresh fixes ? Anyway, it may not be a pleasure process 😨 😵‍💫. I believed that getting pr concentrated is helpful to get merged. People might fear of giant pr, at least, I was hesitated (part of this) to get involved till today.

Could you please take a look at this and alternatives #1848 and #1993 ? @eolivelli @breed @cnauroth @hanm @nkalmar @ztzg @anmolnar @tisonkun @li4wang

@@ -155,11 +155,11 @@ protected void unregisterMetrics() {
}

@Override
public synchronized void shutdown() {
public synchronized void shutdown(boolean fullyShutDown) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a little worried about the modification here has an impact on the invoking chain.

Before modification: Leader.shutdown(String) -> LeaderZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown()
After modification: Leader.shutdown() -> ZooKeeperServer.shutdown()

LeaderZooKeeperServer.shutdown is skipped and containerManager does not stop.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ZooKeeperServer.shutdown() only calls shutdown(false), which is implemented in LeaderZooKeeperServer, and which stops the containerManager. shutdown() isn't overridden anywhere anymore.

@tsuna
Copy link

tsuna commented Jun 13, 2024

Is this PR officially getting superseded by #2152 and #2154?

@jonmv
Copy link
Contributor Author

jonmv commented Jun 13, 2024

Not for me to decide, but I'd think so.

@jonmv
Copy link
Contributor Author

jonmv commented Sep 19, 2024

Superseded by #2111, #2152 and #2154.

@jonmv jonmv closed this Sep 19, 2024
@jonmv jonmv deleted the jonmv/ZOOKEEPER-4541 branch September 19, 2024 07:57
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

10 participants